315c91
@@ -29,6 +29,11 @@
import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -1399,7 +1404,9 @@
implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
   }
 
   /**
-   * We do the following.
+   * We do the following in a different thread.  If it is not completed
+   * in time, we will time it out and assume it is not easy to recover.
+   *
    * 1. Create a new ZK session. (since our current one is expired)
    * 2. Try to become a primary master again
    * 3. Initialize all ZK based system trackers.
@@ -1410,29 +1417,53 @@
implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
    * @return True if we could successfully recover from ZK session expiry.
    * @throws InterruptedException
    * @throws IOException
+   * @throws KeeperException
+   * @throws ExecutionException
    */
   private boolean tryRecoveringExpiredZKSession() throws InterruptedException,
-      IOException, KeeperException {
+      IOException, KeeperException, ExecutionException {
+
     this.zooKeeper = new ZooKeeperWatcher(conf, MASTER + ":"
-        + this.serverName.getPort(), this, true);
+      + this.serverName.getPort(), this, true);
 
-    MonitoredTask status = 
-      TaskMonitor.get().createStatus("Recovering expired ZK session");
-    try {
-      if (!becomeActiveMaster(status)) {
-        return false;
+    Callable<Boolean> callable = new Callable<Boolean> () {
+      public Boolean call() throws InterruptedException,
+          IOException, KeeperException {
+        MonitoredTask status =
+          TaskMonitor.get().createStatus("Recovering expired ZK session");
+        try {
+          if (!becomeActiveMaster(status)) {
+            return Boolean.FALSE;
+          }
+          initializeZKBasedSystemTrackers();
+          // Update in-memory structures to reflect our earlier Root/Meta assignment.
+          assignRootAndMeta(status);
+          // process RIT if any
+          // TODO: Why does this not call AssignmentManager.joinCluster?  Otherwise
+          // we are not processing dead servers if any.
+          assignmentManager.processDeadServersAndRegionsInTransition();
+          return Boolean.TRUE;
+        } finally {
+          status.cleanup();
+        }
+      }
+    };
+
+    long timeout =
+      conf.getLong("hbase.master.zksession.recover.timeout", 300000);
+    java.util.concurrent.ExecutorService executor =
+      Executors.newSingleThreadExecutor();
+    Future<Boolean> result = executor.submit(callable);
+    executor.shutdown();
+    if (executor.awaitTermination(timeout, TimeUnit.MILLISECONDS)
+        && result.isDone()) {
+      Boolean recovered = result.get();
+      if (recovered != null) {
+        return recovered.booleanValue();
       }
-      initializeZKBasedSystemTrackers();
-      // Update in-memory structures to reflect our earlier Root/Meta assignment.
-      assignRootAndMeta(status);
-      // process RIT if any
-      // TODO: Why does this not call AssignmentManager.joinCluster?  Otherwise
-      // we are not processing dead servers if any.
-      this.assignmentManager.processDeadServersAndRegionsInTransition();
-      return true;
-    } finally {
-      status.cleanup();
     }
+    executor.shutdownNow();
+    return false;
   }
 
   /**
